8f273c
@@ -17,24 +17,14 @@
  */
 package org.apache.hadoop.hive.druid.io;
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.metamx.common.lifecycle.Lifecycle;
-import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.HttpClientConfig;
-import com.metamx.http.client.HttpClientInit;
-import io.druid.query.Druids;
-import io.druid.query.Druids.SegmentMetadataQueryBuilder;
-import io.druid.query.Druids.SelectQueryBuilder;
-import io.druid.query.Druids.TimeBoundaryQueryBuilder;
-import io.druid.query.Query;
-import io.druid.query.Result;
-import io.druid.query.metadata.metadata.SegmentAnalysis;
-import io.druid.query.metadata.metadata.SegmentMetadataQuery;
-import io.druid.query.select.PagingSpec;
-import io.druid.query.select.SelectQuery;
-import io.druid.query.spec.MultipleIntervalSegmentSpec;
-import io.druid.query.timeboundary.TimeBoundaryQuery;
-import io.druid.query.timeboundary.TimeBoundaryResultValue;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.calcite.adapter.druid.DruidDateTimeUtils;
 import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.commons.lang3.StringEscapeUtils;
@@ -67,13 +57,28 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.BaseQuery;
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.Druids.SelectQueryBuilder;
+import io.druid.query.Druids.TimeBoundaryQueryBuilder;
+import io.druid.query.Query;
+import io.druid.query.Result;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import io.druid.query.select.PagingSpec;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.spec.MultipleIntervalSegmentSpec;
+import io.druid.query.timeboundary.TimeBoundaryQuery;
+import io.druid.query.timeboundary.TimeBoundaryResultValue;
 
 /**
  * Druid query based input format.
@@ -131,13 +136,19 @@
     JobContext jobContext = ShimLoader.getHadoopShims().newJobContext(job);
     Path[] paths = FileInputFormat.getInputPaths(jobContext);
 
+    // We need to deserialize and serialize query so intervals are written in the JSON
+    // Druid query with user timezone, as this is default Hive time semantics.
+    // Then, create splits with the Druid queries.
     switch (druidQueryType) {
       case Query.TIMESERIES:
       case Query.TOPN:
       case Query.GROUP_BY:
-        return new HiveDruidSplit[] { new HiveDruidSplit(address, druidQuery, paths[0]) };
+        return new HiveDruidSplit[] { new HiveDruidSplit(address,
+                deserializeSerialize(druidQuery), paths[0]) };
       case Query.SELECT:
-        return splitSelectQuery(conf, address, druidQuery, paths[0]);
+        SelectQuery selectQuery = DruidStorageHandlerUtils.JSON_MAPPER.readValue(
+                druidQuery, SelectQuery.class);
+        return splitSelectQuery(conf, address, selectQuery, paths[0]);
       default:
         throw new IOException("Druid query type not recognized");
     }
@@ -158,7 +169,7 @@
private static String createSelectStarQuery(String dataSource) throws IOExceptio
   /* Method that splits Select query depending on the threshold so read can be
    * parallelized */
   private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address,
-          String druidQuery, Path dummyPath
+          SelectQuery query, Path dummyPath
   ) throws IOException {
     final int selectThreshold = (int) HiveConf.getIntVar(
             conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
@@ -166,12 +177,6 @@
private static String createSelectStarQuery(String dataSource) throws IOExceptio
             .getIntVar(conf, HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION);
     final Period readTimeout = new Period(
             HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT));
-    SelectQuery query;
-    try {
-      query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, SelectQuery.class);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
 
     final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false);
     if (isFetch) {
@@ -353,6 +358,13 @@
private static String createSelectStarQuery(String dataSource) throws IOExceptio
     return newIntervals;
   }
 
+  private static String deserializeSerialize(String druidQuery)
+          throws JsonParseException, JsonMappingException, IOException {
+    BaseQuery<?> deserializedQuery = DruidStorageHandlerUtils.JSON_MAPPER.readValue(
+            druidQuery, BaseQuery.class);
+    return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(deserializedQuery);
+  }
+
   @Override
   public org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> getRecordReader(
           org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter
